Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[core][compiled graphs] Overlap computation and communication #47586

Merged
merged 16 commits into from
Oct 31, 2024

Conversation

ruisearch42
Copy link
Contributor

@ruisearch42 ruisearch42 commented Sep 10, 2024

Why are these changes needed?

This PR supports overlapping computation and communication for GPU tasks, as described in https://docs.google.com/document/d/1AkAqrMPadk1rMyjKE4VN4bq058z36fgBcx0i4dHIW20/edit#heading=h.8jw8z0hmgva0

The scope is send/recv but does not include collectives.

Checked perf improvement with test_torch_tensor_dag::test_torch_tensor_nccl_overlap, result is consistent and roughly:
overlap_gpu_communication=False, duration=1.0124679207801819
overlap_gpu_communication=True, duration=0.8186687417328358

Related issue number

Closes #47016
Closes #48191

Visualization

test_torch_tensor_nccl_overlap:

unit_test

zero bubble:

compiled_graph_schedule

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Manual tests with 1f1b and zbh1 schedules, performance is slightly better
    • Release tests
    • This PR is not tested :(

@ruisearch42 ruisearch42 force-pushed the overlap_comm branch 2 times, most recently from 9e13938 to ccb561c Compare September 17, 2024 22:36
@ruisearch42 ruisearch42 changed the title WIP [aDAG] Overlap computation and communication Sep 17, 2024
@ruisearch42 ruisearch42 force-pushed the overlap_comm branch 2 times, most recently from 6b66049 to 4e752b2 Compare September 18, 2024 22:55
@stephanie-wang stephanie-wang self-assigned this Sep 19, 2024
Copy link
Contributor

@stephanie-wang stephanie-wang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Main comment is to try to reuse existing codepaths for executing tasks and reading/writing local args. I think the code will be much more robust this way, plus we need to do it anyway to support enabling overlapping per-task or object.

Seems possible to do if we wrap all inputs/outputs with a wrapper class like this, maybe we need to update the channel reading/writing:

@dataclass
class Value:
  value: Any
  # If set, then reader needs to synchronize before reading value.
  cuda_event: Optional[cp.cuda.Event]

Also should think about how we can unit-test this. Ideally we should try to write a mocked test, maybe something like this one.

python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/context.py Outdated Show resolved Hide resolved
python/ray/experimental/channel/nccl_group.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I really think we need to unify the execution loop. The reason is the test space becomes much larger (we need to also make sure existing case works correctly when overlap is used).

What about we always assume send/recv is unblocking and return Future? And if send/recv is a blocking, the future is returned after wait is finished. Otherwise, it just returns future. It is same as how gloo apis also work

python/ray/dag/context.py Outdated Show resolved Hide resolved
python/ray/dag/constants.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Outdated Show resolved Hide resolved
python/ray/experimental/channel/nccl_group.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/tests/experimental/test_torch_tensor_dag.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

2 additional comments;

  • Before proceeding refactoring, is it possible to unblock @woshiyyya? Maybe one easy solution is to build a custom wheel (you can ping reef team to learn how to do it) so that he can just use it for multi node tests.
  • right now, we have 1 stream per recv/send for 1 process. I wonder if we need a stream per channel or should manage a queue of streams to have multiple nccl channels?

@ruisearch42
Copy link
Contributor Author

  • Before proceeding refactoring, is it possible to unblock @woshiyyya? Maybe one easy solution is to build a custom wheel (you can ping reef team to learn how to do it) so that he can just use it for multi node tests.

Yes that's my plan.

  • right now, we have 1 stream per recv/send for 1 process. I wonder if we need a stream per channel or should manage a queue of streams to have multiple nccl channels?

I think initially we can start with 1 stream. Whether multiple streams can provide better performance would depend on a few factors, and may need a bit of design, although we can perhaps try it.

@rkooo567
Copy link
Contributor

Yeah sounds good. Pp anyway has 1:1 only. We can revisit and benchmark when we have more than 1 input output use cases

@ruisearch42
Copy link
Contributor Author

ruisearch42 commented Sep 20, 2024

Newer commits are for graph visualization prototype, just a draft for now and no need to review new code.

@ruisearch42 ruisearch42 force-pushed the overlap_comm branch 3 times, most recently from a419483 to fc5bbb7 Compare October 4, 2024 22:55
@ruisearch42 ruisearch42 marked this pull request as ready for review October 7, 2024 18:54
@ruisearch42 ruisearch42 added the go add ONLY when ready to merge, run all tests label Oct 7, 2024
Copy link
Contributor

@stephanie-wang stephanie-wang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is better but I believe we can still clean up the GPUFuture interface to make the code more robust. Ideally we want to use .wait() everywhere, not just if isinstance(GPUFuture).

Will take a look at the scheduling and testing code separately.

python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/experimental/channel/gpu_communicator.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Show resolved Hide resolved
python/ray/experimental/channel/nccl_group.py Outdated Show resolved Hide resolved
python/ray/experimental/channel/nccl_group.py Outdated Show resolved Hide resolved
python/ray/experimental/channel/nccl_group.py Show resolved Hide resolved
python/ray/experimental/channel/nccl_group.py Show resolved Hide resolved
python/ray/experimental/channel/shared_memory_channel.py Outdated Show resolved Hide resolved
@kevin85421 kevin85421 self-assigned this Oct 9, 2024
Copy link
Member

@kevin85421 kevin85421 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am still reviewing the PR. All comments with "nit" are not necessary to be addressed in this PR.

python/ray/dag/dag_node_operation.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Show resolved Hide resolved
@ruisearch42 ruisearch42 force-pushed the overlap_comm branch 2 times, most recently from ff439f7 to 00930d2 Compare October 12, 2024 00:42
Copy link
Contributor

@stephanie-wang stephanie-wang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm there might be some misunderstanding about the suggestion to use futures. I wasn't imagining that it would need this much code change, but please let me know if my comments don't make sense.

python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/dag_operation_future.py Outdated Show resolved Hide resolved
python/ray/dag/dag_operation_future.py Outdated Show resolved Hide resolved
python/ray/dag/dag_operation_future.py Outdated Show resolved Hide resolved
python/ray/dag/dag_operation_future.py Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/dag_operation_future.py Outdated Show resolved Hide resolved
@ruisearch42
Copy link
Contributor Author

ruisearch42 commented Oct 12, 2024

Hmm there might be some misunderstanding about the suggestion to use futures. I wasn't imagining that it would need this much code change, but please let me know if my comments don't make sense.

Yeah I think the main challenge is we need to examine write values (exception check, multi-return value extraction) earlier than they are written. Now that these values are futures, we need additional handling: the problematic case is _GPUFuture that needs the send_stream to call wait() with.

Previously I came up with two approaches:

  1. Transformed futures: this is the current approach with WrappedFuture, ListFuture etc.
    This approach wraps the base future and a lambda inside another future, and when the outer future is finally waited on, perform the lambda.

  2. Support access to _NcclGroup._send_stream in earlier phases of the data flow (e.g., in the execution loop operations rather than nccl_group.send()), so that we can make the _send_stream wait on the future earlier
    I think this is doable if e.g., in ExecutableTask we can access the _send_stream that is to be used in send(). And I figured out a code path, the logic is roughly:

  • nccl_group_id created when the nccl group is created at compile time
  • Nccl_group_id is set to each TorchTensorType, which is further populated to TorchTensorNcclChannel
  • nccl_group_id is used to retrieve the _NcclGroup from current_context()
  • With _NcclGroup we can access _send_stream
  • nccl_group_id can be passed via this path: ExecutableTask <- CompiledTask <- DAGNode <- _type_hint

I think approach2 leaves out changes to the channel interface but leaks the NcclGroup send_stream implementation details, and approach1 touches a lot of interfaces but things are consistent. Originally I thought about implementing approach2 but found approach1 may be a bit cleaner, although it changes the interfaces quite a bit.

Let me know how you think about it.

@stephanie-wang
Copy link
Contributor

Now that these values are futures, we need additional handling: the problematic case is _GPUFuture that needs the send_stream to call wait() with.

This can be fixed by passing send_stream through the context. I saw your comment about _send_stream not being the current stream but what I meant is that we should change other parts of the code to make sure it is the current stream. I.e. right before this line, set current stream to _send_stream, _recv_stream, etc.

2. Support access to `_NcclGroup._send_stream` in earlier phases of the data flow (e.g., in the execution loop operations rather than nccl_group.send()), so that we can make the `_send_stream` wait on the future earlier

I don't understand why we need to do the wait earlier. It should be okay to just do the wait right before starting to execute the _DAGNodeOperation, same as right now. The stream wait on the event is asynchronous so it shouldn't be a performance issue.

@ruisearch42
Copy link
Contributor Author

I don't understand why we need to do the wait earlier. It should be okay to just do the wait right before starting to execute the _DAGNodeOperation, same as right now. The stream wait on the event is asynchronous so it shouldn't be a performance issue.

Yeah what I meant by "earlier" in

Support access to `_NcclGroup._send_stream` in earlier phases of the data flow (e.g., in the execution loop operations rather than nccl_group.send()), so that we can make the `_send_stream` wait on the future earlier

is exactly this line, and it needs the _send_stream to be passed via ExecutableTask, roughly with this logic I mentioned in approach2:

nccl_group_id created when the nccl group is created at compile time
nccl_group_id is set to each TorchTensorType, which is further populated to TorchTensorNcclChannel
nccl_group_id is used to retrieve the _NcclGroup from current_context()
With _NcclGroup we can access _send_stream
nccl_group_id can be passed via this path: ExecutableTask <- CompiledTask <- DAGNode <- _type_hint

So I think approach2 is what you preferred? If so, I can change to implement it.

@stephanie-wang
Copy link
Contributor

I don't understand why we need to do the wait earlier. It should be okay to just do the wait right before starting to execute the _DAGNodeOperation, same as right now. The stream wait on the event is asynchronous so it shouldn't be a performance issue.

Yeah what I meant by "earlier" in

Support access to `_NcclGroup._send_stream` in earlier phases of the data flow (e.g., in the execution loop operations rather than nccl_group.send()), so that we can make the `_send_stream` wait on the future earlier

is exactly this line, and it needs the _send_stream to be passed via ExecutableTask, roughly with this logic I mentioned in approach2:

nccl_group_id created when the nccl group is created at compile time
nccl_group_id is set to each TorchTensorType, which is further populated to TorchTensorNcclChannel
nccl_group_id is used to retrieve the _NcclGroup from current_context()
With _NcclGroup we can access _send_stream
nccl_group_id can be passed via this path: ExecutableTask <- CompiledTask <- DAGNode <- _type_hint

So I think approach2 is what you preferred? If so, I can change to implement it.

Ah okay, got it, thanks for the explanation. Yes just to be clear, I prefer approach2 for the following reasons:

  1. we don't leak details of overlapping communication etc outside of the DAGNodeOperation
  2. we can call .wait() in only one place (better for testing/robustness)
  3. simpler interfaces; i.e. we only need GPUFuture vs a future that's already ready, and don't need the WrappedFuture, ListFuture, etc.

Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
ruisearch42 and others added 2 commits October 27, 2024 17:49
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
up
Signed-off-by: Rui Qiao <ubuntu@ip-172-31-15-128.us-west-2.compute.internal>
Rui Qiao and others added 5 commits October 29, 2024 17:35
up
Signed-off-by: Rui Qiao <ubuntu@ip-172-31-15-128.us-west-2.compute.internal>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
up
Signed-off-by: Rui Qiao <ubuntu@ip-172-31-15-128.us-west-2.compute.internal>
up
Signed-off-by: Rui Qiao <ubuntu@ip-172-31-15-128.us-west-2.compute.internal>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
)


class AbstractNcclGroup(GPUCommunicator):
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Need to move this from test_collective_dag.py, otherwise there is the following error:

>                       raise value.as_instanceof_cause()
E                       ray.exceptions.RayTaskError(RaySystemError): ray::CPUTorchTensorWorker.__ray_call__() (pid=2553478, ip=172.31.15.128, actor_id=a378363d53bded77aa33e28301000000, repr=<test_collective_dag.CPUTorchTensorWorker object at 0x7f02403f7c40>)
E                         At least one of the input arguments for this task could not be computed:
E                       ray.exceptions.RaySystemError: System error: No module named 'test_collective_dag'
E                       traceback: Traceback (most recent call last):
E                       ModuleNotFoundError: No module named 'test_collective_dag'

Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. One last request for unit tests

python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/dag_operation_future.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node_operation.py Show resolved Hide resolved
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
Signed-off-by: Rui Qiao <ruisearch42@gmail.com>
@rkooo567
Copy link
Contributor

lmk when ti is ready to merge! super excited for this feature.

After this can you follow up

  • design for cross dag overlap
  • release tests for overlap compute/comm

@ruisearch42 ruisearch42 changed the title [aDAG] Overlap computation and communication [core][compiled graphs] Overlap computation and communication Oct 31, 2024
@rkooo567 rkooo567 merged commit 0d94d13 into ray-project:master Oct 31, 2024
5 checks passed
Jay-ju pushed a commit to Jay-ju/ray that referenced this pull request Nov 5, 2024
…oject#47586)

This PR supports overlapping computation and communication for GPU tasks, as described in https://docs.google.com/document/d/1AkAqrMPadk1rMyjKE4VN4bq058z36fgBcx0i4dHIW20/edit#heading=h.8jw8z0hmgva0

The scope is send/recv but does not include collectives.

Checked perf improvement with test_torch_tensor_dag::test_torch_tensor_nccl_overlap, result is consistent and roughly:
overlap_gpu_communication=False, duration=1.0124679207801819
overlap_gpu_communication=True, duration=0.8186687417328358
JP-sDEV pushed a commit to JP-sDEV/ray that referenced this pull request Nov 14, 2024
…oject#47586)

This PR supports overlapping computation and communication for GPU tasks, as described in https://docs.google.com/document/d/1AkAqrMPadk1rMyjKE4VN4bq058z36fgBcx0i4dHIW20/edit#heading=h.8jw8z0hmgva0

The scope is send/recv but does not include collectives.

Checked perf improvement with test_torch_tensor_dag::test_torch_tensor_nccl_overlap, result is consistent and roughly:
overlap_gpu_communication=False, duration=1.0124679207801819
overlap_gpu_communication=True, duration=0.8186687417328358
Signed-off-by: JP-sDEV <jon.pablo80@gmail.com>
mohitjain2504 pushed a commit to mohitjain2504/ray that referenced this pull request Nov 15, 2024
…oject#47586)

This PR supports overlapping computation and communication for GPU tasks, as described in https://docs.google.com/document/d/1AkAqrMPadk1rMyjKE4VN4bq058z36fgBcx0i4dHIW20/edit#heading=h.8jw8z0hmgva0

The scope is send/recv but does not include collectives.

Checked perf improvement with test_torch_tensor_dag::test_torch_tensor_nccl_overlap, result is consistent and roughly:
overlap_gpu_communication=False, duration=1.0124679207801819
overlap_gpu_communication=True, duration=0.8186687417328358

Signed-off-by: mohitjain2504 <mohit.jain@dream11.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
4 participants